KAFKA-20115: Group coordinator fails to unload metadata when no longer leader or follower#21396
KAFKA-20115: Group coordinator fails to unload metadata when no longer leader or follower#21396brandboat wants to merge 4 commits intoapache:4.2from
Conversation
…r leader or follower Signed-off-by: Kuan-Po Tseng <brandboat@gmail.com>
|
cc @clolov This is a potential blocker for 4.2. |
I gave him the heads-up about the bad news offline. Hopefully, he's still my fried after this. 😢 |
squah-confluent
left a comment
There was a problem hiding this comment.
Thank you for the fix!
Some initial thoughts:
| } | ||
|
|
||
| @Test | ||
| public void testScheduleUnloadingWithPendingBatchWhenPartitionWriterConfigThrows() { |
There was a problem hiding this comment.
Could we move this together with the rest of the testScheduleUnloading* tests?
| new CoordinatorRuntime.Builder<MockCoordinatorShard, String>() | ||
| .withTime(timer.time()) | ||
| .withTimer(timer) | ||
| .withDefaultWriteTimeOut(Duration.ofMillis(20)) |
There was a problem hiding this comment.
nit: Could we use DEFAULT_WRITE_TIMEOUT here unless there's a reason to use a different timeout?
| .withDefaultWriteTimeOut(Duration.ofMillis(20)) | |
| .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) |
Signed-off-by: Kuan-Po Tseng <brandboat@gmail.com>
| } | ||
|
|
||
| @Test | ||
| public void testScheduleUnloadingWithPendingBatchWhenPartitionWriterConfigThrows() { |
There was a problem hiding this comment.
I'm thinking about adding integration tests. For example:
@ClusterTest(
brokers = 2,
types = {Type.KRAFT},
serverProperties = {
@ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
@ClusterConfigProperty(key = "group.coordinator.append.linger.ms", value = "3000")
}
)
public void test(ClusterInstance clusterInstance) throws InterruptedException, ExecutionException, TimeoutException {
try (var producer = clusterInstance.<byte[], byte[]>producer()) {
producer.send(new ProducerRecord<>("topic", "value".getBytes(StandardCharsets.UTF_8)));
}
try (var admin = clusterInstance.admin()) {
admin.createTopics(List.of(new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, Map.of(0, List.of(0))))).all().get();
}
try (var consumer = clusterInstance.consumer(Map.of(ConsumerConfig.GROUP_ID_CONFIG, "test-group"));
var admin = clusterInstance.admin()) {
consumer.subscribe(List.of("topic"));
while (consumer.poll(Duration.ofMillis(100)).isEmpty()) {
// empty body
}
// append records to coordinator
consumer.commitSync();
// unload the coordinator by changing leader (0 -> 1)
admin.alterPartitionReassignments(Map.of(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0),
Optional.of(new NewPartitionReassignment(List.of(1))))).all().get();
}
Function<GroupCoordinator, List<TopicPartition>> partitionsInGroupMetrics = service -> assertDoesNotThrow(() -> {
var f0 = GroupCoordinatorService.class.getDeclaredField("groupCoordinatorMetrics");
f0.setAccessible(true);
var f1 = GroupCoordinatorMetrics.class.getDeclaredField("shards");
f1.setAccessible(true);
return List.copyOf(((Map<TopicPartition, ?>) f1.get(f0.get(service))).keySet());
});
// the offset partition should NOT be hosted by multiple coordinators
var tps = clusterInstance.brokers().values().stream()
.flatMap(b -> partitionsInGroupMetrics.apply(b.groupCoordinator()).stream()).toList();
assertEquals(1, tps.size());
}WDYT?
There was a problem hiding this comment.
Sure, why not? Thanks for the thorough integration test!
Signed-off-by: Kuan-Po Tseng <brandboat@gmail.com>
| deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception()); | ||
| failCurrentBatch(Errors.NOT_COORDINATOR.exception()); | ||
| // There is no need to free the current batch, as we will be closing all related resources anyway. | ||
| failCurrentBatch(Errors.NOT_COORDINATOR.exception(), false); |
There was a problem hiding this comment.
Would you mind adding a readable method? for example:
private void failCurrentBatchWithoutRelease(Throwable t) {
failCurrentBatch(t, false);
}| @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, value = "3000") | ||
| } | ||
| ) | ||
| public void testSingleCoordinatorOwnershipAfterPartitionReassignment(ClusterInstance clusterInstance) throws InterruptedException, ExecutionException, TimeoutException { |
There was a problem hiding this comment.
@brandboat could you open a PR against trunk to improve the test coverage?
Signed-off-by: Kuan-Po Tseng <brandboat@gmail.com>
When a broker loses leadership of a __consumer_offsets partition while a
write batch is pending, the coordinator unload process fails because
freeCurrentBatch() attempts to access partition writer configuration
which throws NOT_LEADER_OR_FOLLOWER exception.
This commit fixes the issue by skipping buffer release during unload
since all related resources are being closed anyway.